Skip to content

[fix][broker] Correct two race conditions in the tracker code and logic bug in InMemoryDelayedDeliveryTracker that failed with NoSuchElementException#25681

Merged
lhotari merged 7 commits into
apache:masterfrom
chamons:broker_race_condition_fix
May 8, 2026

Conversation

@chamons
Copy link
Copy Markdown
Contributor

@chamons chamons commented May 5, 2026

Fixes #25617

Motivation

While running a production system under load, we found a traffic pattern that could cause uncaught exceptions in the brokers of our pulsar cluster. If you have a large spike of messages with a matching deliver_at_time, it is possible to trigger java.util.NoSuchElementException and other exceptions:

	at it.unimi.dsi.fastutil.longs.Long2ObjectAVLTreeMap.firstLongKey(Long2ObjectAVLTreeMap.java:934) ~[it.unimi.dsi-fastutil-8.5.16.jar:?]
	at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.nextDeliveryTime(InMemoryDelayedDeliveryTracker.java:273) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2]
	at org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker.updateTimer(AbstractDelayedDeliveryTracker.java:99) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2]
	at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.getScheduledMessages(InMemoryDelayedDeliveryTracker.java:229) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2]
	at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.getMessagesToReplayNow(PersistentDispatcherMultipleConsumers.java:1327) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2]
	at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readMoreEntries(PersistentDispatcherMultipleConsumers.java:385) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2]
	at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.lambda$readMoreEntriesAsync$4(PersistentDispatcherMultipleConsumers.java:324) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2]

This is a significant issue, as exceptions here were uncaught and put the broker into an invalid state that prevented some delayed messages from being delivered until restart.

Using our external repro as a guide, we were able to isolate two race conditions and a logic bug that triggered exceptions that matched these.

With these three fixes, we were able to run our sample case for 30 minutes straight without exceptions (before hand it would trigger within 3-5 minutes).

Modifications

Posting each commit's description here for simplicity:

[fix][broker] Correct multiple race conditions in PersistentDispatcherMultipleConsumers
- https://github.com/apache/pulsar/issues/25617

DelayedDeliveryTracker is not thread safe, and any access to it must be done
holding the object lock. There were five cases I found, four of them I could
correct just by adding synchronized to the method declaration. In one of them
I used a manual scope since it was only a subset of the method.

This is a significant issue, as exceptions here were uncaught and put
the broker into an invalid state that prevented some delayed messages
from being delivered until restart.

The included unit test failed 100% of the time when run locally without the fix.
[fix][broker] Fix multiple race conditions in InMemoryDelayedDeliveryTracker

While reviewing the code for more issues to explain the crashes we were still seeing,
I noticed that InMemoryDelayedDeliveryTracker contains two major state fields:

- delayedMessageMap
- delayedMessagesCount

These are kept in sync in various methods, but those methods were _not_ correctly
synchronized, which meant it was trivial for us to get them out of sync
and then crash peeking at the map.

Added a unit test that triggers 100% of the time before the fix.
[fix][broker] Fix crash when duplicate ledger/entry messages come into InMemoryDelayedDeliveryTracker

When the same ledger/entry messages come into InMemoryDelayedDeliveryTracker, we can desync the
delayedMessagesCount/delayedMessageMap count such that there are less items in the map than the count

Instead, we track if the bitmap doesn't already contain the item, and only increment delayedMessagesCount if not.

Verifying this change

  • Make sure that the change passes the CI checks.

I have a clean CI here

image

The included unit tests fail 100% of the time when run locally without the fix, and we were able to run https://github.com/chamons/pulsar-scheduled-exception-repro without issues with the fixes included.

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment

@chamons
Copy link
Copy Markdown
Contributor Author

chamons commented May 6, 2026

@merlimat - We found that this is only a partial fix for #25617, as there is at least one other set of race conditions that can trigger it.

Would you prefer a single larger PR with all of the included or should I fix this up and land piecemeal?

@chamons chamons force-pushed the broker_race_condition_fix branch 3 times, most recently from 39814a6 to c11c32f Compare May 7, 2026 19:14
@chamons chamons changed the title [fix][broker] Correct multiple race conditions in PersistentDispatcherMultipleConsumers [fix][broker] Correct two race conditions in the tracker code and logic bug in InMemoryDelayedDeliveryTracker that failed with NoSuchElementException May 7, 2026
@chamons chamons force-pushed the broker_race_condition_fix branch 2 times, most recently from 992ab28 to 0038cf4 Compare May 7, 2026 19:42
chamons added 3 commits May 7, 2026 15:19
…rMultipleConsumers

- apache#25617

DelayedDeliveryTracker is not thread safe, and any access to it must be done
holding the object lock. There were five cases I found, four of them I could
correct just by adding synchronized to the method declaration. In one of them
I used a manual scope since it was only a subset of the method.

This is a significant issue, as exceptions here were uncaught and put
the broker into an invalid state that prevented some delayed messages
from being delivered until restart.

The included unit test failed 100% of the time when run locally without the fix.
…Tracker

While reviewing the code for more issues to explain the crashes we were still seeing,
I noticed that InMemoryDelayedDeliveryTracker contains two major state fields:

- delayedMessageMap
- delayedMessagesCount

These are kept in sync in various methods, but those methods were _not_ correctly
synchronized, which meant it was trivial for us to get them out of sync
and then crash peeking at the map.

Added a unit test that triggers 100% of the time before the fix.
…o InMemoryDelayedDeliveryTracker

When the same ledger/entry messages come into InMemoryDelayedDeliveryTracker, we can desync the
delayedMessagesCount/delayedMessageMap count such that there are less items in the map than the count

If then we pop down to zero (not clear), we can get:

```
java.util.NoSuchElementException
        at java.base/java.util.TreeMap.key(TreeMap.java:1637)
        at java.base/java.util.TreeMap.firstKey(TreeMap.java:302)
        at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.nextDeliveryTime(InMemoryDelayedDeliveryTracker.java:291)
        at org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker.updateTimer(AbstractDelayedDeliveryTracker.java:117)
        at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.getScheduledMessages(InMemoryDelayedDeliveryTracker.java:247)
        at org.apache.pulsar.broker.delayed.InMemoryDeliveryTrackerTest.testAddMultipleMessagesSameWindow(InMemoryDeliveryTrackerTest.java:328)
```

Instead, we track if the bitmap doesn't already contain the item, and only increment delayedMessagesCount if not.
@chamons chamons force-pushed the broker_race_condition_fix branch from 0038cf4 to 66df8a9 Compare May 7, 2026 20:19
@chamons chamons marked this pull request as ready for review May 7, 2026 21:12
@lhotari lhotari added this to the 5.0.0-M1 milestone May 8, 2026
@chamons chamons requested review from lhotari and merlimat May 8, 2026 16:46
@chamons
Copy link
Copy Markdown
Contributor Author

chamons commented May 8, 2026

I have resolved all current PR review comments and am ready for a re-review.

Copy link
Copy Markdown
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks for the important bug fix and contribution @chamons! Good job!

@lhotari lhotari merged commit 6cbf4d2 into apache:master May 8, 2026
43 checks passed
@chamons
Copy link
Copy Markdown
Contributor Author

chamons commented May 8, 2026

Thanks @lhotari - Do we have any idea when this fix might make it into a public release? Could it make it into a 4.x branch?

We'd like to make some plans, and knowing what timetable to expect would be really helpful.

@merlimat
Copy link
Copy Markdown
Contributor

merlimat commented May 8, 2026

Yes, it's a bug fix and it will be backported into 4.0.x (prob 4.0.11) release

lhotari pushed a commit that referenced this pull request May 11, 2026
…ic bug in InMemoryDelayedDeliveryTracker that failed with NoSuchElementException (#25681)

(cherry picked from commit 6cbf4d2)
lhotari pushed a commit that referenced this pull request May 11, 2026
…ic bug in InMemoryDelayedDeliveryTracker that failed with NoSuchElementException (#25681)

(cherry picked from commit 6cbf4d2)
lhotari pushed a commit that referenced this pull request May 11, 2026
…ic bug in InMemoryDelayedDeliveryTracker that failed with NoSuchElementException (#25681)

(cherry picked from commit 6cbf4d2)
@lhotari
Copy link
Copy Markdown
Member

lhotari commented May 11, 2026

@chamons I have cherry-picked in this to branch-4.0, branch-4.1 and branch-4.2 . Since the implementations in each branch is slightly different, the change has been backported. Please note that 4.1.x is no longer supported, so the release won't be made on that branch although the fix is present on the branch. It's recommended to upgrade to 4.2.2 once it comes out within about a month. In the mean time, you can build your own release. The Pulsar release is performed using these steps: https://pulsar.apache.org/contribute/release-process/#build-release-artifacts . The "Build release artifacts" and "Stage docker images" are the only required steps in this case.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug] NoSuchElementException in InMemoryDelayedDeliveryTracker.nextDeliveryTime broke message dispatching until broker was restarted

3 participants